/
scopeRemoteRunTypes.go
559 lines (499 loc) · 16 KB
/
scopeRemoteRunTypes.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
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
// Copyright 2021 Matrix Origin
//
// 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package compile
import (
"context"
"fmt"
"hash/crc32"
"runtime"
"sync/atomic"
"time"
"github.com/matrixorigin/matrixone/pkg/logservice"
"github.com/google/uuid"
"github.com/matrixorigin/matrixone/pkg/cnservice/cnclient"
"github.com/matrixorigin/matrixone/pkg/common/moerr"
"github.com/matrixorigin/matrixone/pkg/common/morpc"
"github.com/matrixorigin/matrixone/pkg/common/mpool"
"github.com/matrixorigin/matrixone/pkg/container/batch"
"github.com/matrixorigin/matrixone/pkg/container/types"
"github.com/matrixorigin/matrixone/pkg/defines"
"github.com/matrixorigin/matrixone/pkg/fileservice"
"github.com/matrixorigin/matrixone/pkg/lockservice"
"github.com/matrixorigin/matrixone/pkg/logutil"
"github.com/matrixorigin/matrixone/pkg/pb/pipeline"
"github.com/matrixorigin/matrixone/pkg/pb/plan"
"github.com/matrixorigin/matrixone/pkg/perfcounter"
"github.com/matrixorigin/matrixone/pkg/queryservice"
"github.com/matrixorigin/matrixone/pkg/sql/colexec"
"github.com/matrixorigin/matrixone/pkg/txn/client"
"github.com/matrixorigin/matrixone/pkg/udf"
"github.com/matrixorigin/matrixone/pkg/vm/engine"
"github.com/matrixorigin/matrixone/pkg/vm/process"
)
const (
maxMessageSizeToMoRpc = 64 * mpool.MB
// HandleNotifyTimeout
// todo: this is a bad design here.
// we should do the waiting work in the prepare stage of the dispatch operator but not in the exec stage.
// do the waiting work in the exec stage can save some execution time, but it will cause an unstable waiting time.
// (because we cannot control the execution time of the running sql,
// and the coming time of the first batch of the result is not a constant time.)
// see the codes in pkg/sql/colexec/dispatch/dispatch.go:waitRemoteRegsReady()
//
// need to fix this in the future. for now, increase it to make tpch1T can run on 3 CN
HandleNotifyTimeout = 300 * time.Second
)
// cnInformation records service information to help handle messages.
type cnInformation struct {
cnAddr string
storeEngine engine.Engine
fileService fileservice.FileService
lockService lockservice.LockService
queryService queryservice.QueryService
hakeeper logservice.CNHAKeeperClient
udfService udf.Service
aicm *defines.AutoIncrCacheManager
}
// processHelper records source process information to help
// rebuild the process at the remote node.
type processHelper struct {
id string
lim process.Limitation
unixTime int64
accountId uint32
txnOperator client.TxnOperator
txnClient client.TxnClient
sessionInfo process.SessionInfo
analysisNodeList []int32
}
// messageSenderOnClient is a structure
// for sending message and receiving results on cn-client.
type messageSenderOnClient struct {
ctx context.Context
ctxCancel context.CancelFunc
streamSender morpc.Stream
receiveCh chan morpc.Message
c *Compile
}
func newMessageSenderOnClient(
ctx context.Context, c *Compile, toAddr string) (*messageSenderOnClient, error) {
var sender = new(messageSenderOnClient)
streamSender, err := cnclient.GetStreamSender(toAddr)
if err != nil {
return sender, err
}
sender.streamSender = streamSender
if _, ok := ctx.Deadline(); !ok {
sender.ctx, sender.ctxCancel = context.WithTimeout(ctx, time.Second*10000)
} else {
sender.ctx = ctx
}
sender.c = c
return sender, nil
}
// XXX we can set a scope as argument directly next day.
func (sender *messageSenderOnClient) send(
scopeData, procData []byte, messageType pipeline.Method) error {
sdLen := len(scopeData)
if sdLen <= maxMessageSizeToMoRpc {
message := cnclient.AcquireMessage()
message.SetID(sender.streamSender.ID())
message.SetMessageType(pipeline.Method_PipelineMessage)
message.SetData(scopeData)
message.SetProcData(procData)
message.SetSequence(0)
message.SetSid(pipeline.Status_Last)
return sender.streamSender.Send(sender.ctx, message)
}
start := 0
cnt := uint64(0)
for start < sdLen {
end := start + maxMessageSizeToMoRpc
message := cnclient.AcquireMessage()
message.SetID(sender.streamSender.ID())
message.SetMessageType(pipeline.Method_PipelineMessage)
message.SetSequence(cnt)
if end >= sdLen {
message.SetData(scopeData[start:sdLen])
message.SetProcData(procData)
message.SetSid(pipeline.Status_Last)
} else {
message.SetData(scopeData[start:end])
message.SetSid(pipeline.Status_WaitingNext)
}
if err := sender.streamSender.Send(sender.ctx, message); err != nil {
return err
}
cnt++
start = end
}
return nil
}
func (sender *messageSenderOnClient) receiveMessage() (morpc.Message, error) {
select {
case <-sender.ctx.Done():
return nil, nil
case val, ok := <-sender.receiveCh:
if !ok || val == nil {
// ch close
return nil, moerr.NewStreamClosed(sender.ctx)
}
return val, nil
}
}
func (sender *messageSenderOnClient) receiveBatch() (bat *batch.Batch, over bool, err error) {
var val morpc.Message
var m *pipeline.Message
var dataBuffer []byte
for {
val, err = sender.receiveMessage()
if err != nil {
return nil, false, err
}
if val == nil {
return nil, true, nil
}
m = val.(*pipeline.Message)
if info, get := m.TryToGetMoErr(); get {
return nil, false, info
}
if m.IsEndMessage() {
anaData := m.GetAnalyse()
if len(anaData) > 0 {
ana := new(pipeline.AnalysisList)
if err = ana.Unmarshal(anaData); err != nil {
return nil, false, err
}
mergeAnalyseInfo(sender.c.anal, ana)
}
return nil, true, nil
}
if dataBuffer == nil {
dataBuffer = m.Data
} else {
dataBuffer = append(dataBuffer, m.Data...)
}
if m.WaitingNextToMerge() {
continue
}
if m.Checksum != crc32.ChecksumIEEE(dataBuffer) {
return nil, false, moerr.NewInternalErrorNoCtx("Packages delivered by morpc is broken")
}
bat, err = decodeBatch(sender.c.proc.Mp(), sender.c.proc, dataBuffer)
if err != nil {
return nil, false, err
}
return bat, false, nil
}
}
func mergeAnalyseInfo(target *anaylze, ana *pipeline.AnalysisList) {
source := ana.List
if len(target.analInfos) != len(source) {
return
}
for i := range target.analInfos {
n := source[i]
atomic.AddInt64(&target.analInfos[i].OutputSize, n.OutputSize)
atomic.AddInt64(&target.analInfos[i].OutputRows, n.OutputRows)
atomic.AddInt64(&target.analInfos[i].InputRows, n.InputRows)
atomic.AddInt64(&target.analInfos[i].InputSize, n.InputSize)
atomic.AddInt64(&target.analInfos[i].MemorySize, n.MemorySize)
target.analInfos[i].MergeArray(n)
atomic.AddInt64(&target.analInfos[i].TimeConsumed, n.TimeConsumed)
atomic.AddInt64(&target.analInfos[i].WaitTimeConsumed, n.WaitTimeConsumed)
atomic.AddInt64(&target.analInfos[i].DiskIO, n.DiskIO)
atomic.AddInt64(&target.analInfos[i].S3IOByte, n.S3IOByte)
atomic.AddInt64(&target.analInfos[i].S3IOInputCount, n.S3IOInputCount)
atomic.AddInt64(&target.analInfos[i].S3IOOutputCount, n.S3IOOutputCount)
atomic.AddInt64(&target.analInfos[i].NetworkIO, n.NetworkIO)
atomic.AddInt64(&target.analInfos[i].ScanTime, n.ScanTime)
atomic.AddInt64(&target.analInfos[i].InsertTime, n.InsertTime)
}
}
func (sender *messageSenderOnClient) close() {
if sender.ctxCancel != nil {
sender.ctxCancel()
}
// XXX not a good way to deal it if close failed.
_ = sender.streamSender.Close(true)
}
// messageReceiverOnServer is a structure
// for processing received message and writing results back at cn-server.
type messageReceiverOnServer struct {
ctx context.Context
messageId uint64
messageTyp pipeline.Method
messageUuid uuid.UUID
cnInformation cnInformation
// information to build a process.
procBuildHelper processHelper
clientSession morpc.ClientSession
messageAcquirer func() morpc.Message
maxMessageSize int
scopeData []byte
// XXX what's that. So confused.
sequence uint64
// result.
finalAnalysisInfo []*process.AnalyzeInfo
}
func newMessageReceiverOnServer(
ctx context.Context,
cnAddr string,
m *pipeline.Message,
cs morpc.ClientSession,
messageAcquirer func() morpc.Message,
storeEngine engine.Engine,
fileService fileservice.FileService,
lockService lockservice.LockService,
queryService queryservice.QueryService,
hakeeper logservice.CNHAKeeperClient,
udfService udf.Service,
txnClient client.TxnClient,
aicm *defines.AutoIncrCacheManager) messageReceiverOnServer {
receiver := messageReceiverOnServer{
ctx: ctx,
messageId: m.GetId(),
messageTyp: m.GetCmd(),
clientSession: cs,
messageAcquirer: messageAcquirer,
maxMessageSize: maxMessageSizeToMoRpc,
sequence: 0,
}
receiver.cnInformation = cnInformation{
cnAddr: cnAddr,
storeEngine: storeEngine,
fileService: fileService,
lockService: lockService,
queryService: queryService,
hakeeper: hakeeper,
udfService: udfService,
aicm: aicm,
}
switch m.GetCmd() {
case pipeline.Method_PrepareDoneNotifyMessage:
opUuid, err := uuid.FromBytes(m.GetUuid())
if err != nil {
logutil.Errorf("decode uuid from pipeline.Message failed, bytes are %v", m.GetUuid())
panic("cn receive a message with wrong uuid bytes")
}
receiver.messageUuid = opUuid
case pipeline.Method_PipelineMessage:
var err error
receiver.procBuildHelper, err = generateProcessHelper(m.GetProcInfoData(), txnClient)
if err != nil {
logutil.Errorf("decode process info from pipeline.Message failed, bytes are %v", m.GetProcInfoData())
panic("cn receive a message with wrong process bytes")
}
receiver.scopeData = m.Data
default:
logutil.Errorf("unknown cmd %d for pipeline.Message", m.GetCmd())
panic("unknown message type")
}
return receiver
}
func (receiver *messageReceiverOnServer) acquireMessage() (*pipeline.Message, error) {
message, ok := receiver.messageAcquirer().(*pipeline.Message)
if !ok {
return nil, moerr.NewInternalError(receiver.ctx, "get a message with wrong type.")
}
message.SetID(receiver.messageId)
return message, nil
}
// newCompile make and return a new compile to run a pipeline.
func (receiver *messageReceiverOnServer) newCompile() *Compile {
// compile is almost surely wanting a small or middle pool. Later.
mp, err := mpool.NewMPool("compile", 0, mpool.NoFixed)
if err != nil {
panic(err)
}
pHelper, cnInfo := receiver.procBuildHelper, receiver.cnInformation
proc := process.New(
receiver.ctx,
mp,
pHelper.txnClient,
pHelper.txnOperator,
cnInfo.fileService,
cnInfo.lockService,
cnInfo.queryService,
cnInfo.hakeeper,
cnInfo.udfService,
cnInfo.aicm)
proc.UnixTime = pHelper.unixTime
proc.Id = pHelper.id
proc.Lim = pHelper.lim
proc.SessionInfo = pHelper.sessionInfo
proc.SessionInfo.StorageEngine = cnInfo.storeEngine
proc.AnalInfos = make([]*process.AnalyzeInfo, len(pHelper.analysisNodeList))
for i := range proc.AnalInfos {
proc.AnalInfos[i] = process.NewAnalyzeInfo()
proc.AnalInfos[i].NodeId = pHelper.analysisNodeList[i]
}
proc.DispatchNotifyCh = make(chan process.WrapCs)
c := &Compile{
proc: proc,
e: cnInfo.storeEngine,
anal: &anaylze{analInfos: proc.AnalInfos},
addr: receiver.cnInformation.cnAddr,
}
c.proc.Ctx = perfcounter.WithCounterSet(c.proc.Ctx, &c.counterSet)
c.ctx = defines.AttachAccountId(c.proc.Ctx, pHelper.accountId)
c.fill = func(_ any, b *batch.Batch) error {
return receiver.sendBatch(b)
}
c.runtimeFilterReceiverMap = make(map[int32]*runtimeFilterReceiver)
return c
}
func (receiver *messageReceiverOnServer) sendError(
errInfo error) error {
message, err := receiver.acquireMessage()
if err != nil {
return err
}
message.SetID(receiver.messageId)
message.SetSid(pipeline.Status_MessageEnd)
if errInfo != nil {
message.SetMoError(receiver.ctx, errInfo)
}
return receiver.clientSession.Write(receiver.ctx, message)
}
func (receiver *messageReceiverOnServer) sendBatch(
b *batch.Batch) error {
// there's no need to send the nil batch.
if b == nil {
return nil
}
// There is still a memory problem here. If row count is very small, but the cap of batch's vectors is very large,
// to encode will allocate a large memory.
// but I'm not sure how string type store data in vector, so I can't do a simple optimization like vec.col = vec.col[:len].
data, err := types.Encode(b)
if err != nil {
return err
}
checksum := crc32.ChecksumIEEE(data)
dataLen := len(data)
if dataLen <= receiver.maxMessageSize {
m, errA := receiver.acquireMessage()
if errA != nil {
return errA
}
m.SetMessageType(pipeline.Method_BatchMessage)
m.SetData(data)
// XXX too bad.
m.SetCheckSum(checksum)
m.SetSequence(receiver.sequence)
m.SetSid(pipeline.Status_Last)
receiver.sequence++
return receiver.clientSession.Write(receiver.ctx, m)
}
// if data is too large, cut and send
for start, end := 0, 0; start < dataLen; start = end {
m, errA := receiver.acquireMessage()
if errA != nil {
return errA
}
end = start + receiver.maxMessageSize
if end >= dataLen {
end = dataLen
m.SetSid(pipeline.Status_Last)
m.SetCheckSum(checksum)
} else {
m.SetSid(pipeline.Status_WaitingNext)
}
m.SetMessageType(pipeline.Method_BatchMessage)
m.SetData(data[start:end])
m.SetSequence(receiver.sequence)
receiver.sequence++
if errW := receiver.clientSession.Write(receiver.ctx, m); errW != nil {
return errW
}
}
return nil
}
func (receiver *messageReceiverOnServer) sendEndMessage() error {
message, err := receiver.acquireMessage()
if err != nil {
return err
}
message.SetSid(pipeline.Status_MessageEnd)
message.SetID(receiver.messageId)
message.SetMessageType(receiver.messageTyp)
analysisInfo := receiver.finalAnalysisInfo
if len(analysisInfo) > 0 {
anas := &pipeline.AnalysisList{
List: make([]*plan.AnalyzeInfo, len(analysisInfo)),
}
for i, a := range analysisInfo {
anas.List[i] = convertToPlanAnalyzeInfo(a)
}
data, err := anas.Marshal()
if err != nil {
return err
}
message.SetAnalysis(data)
}
return receiver.clientSession.Write(receiver.ctx, message)
}
func generateProcessHelper(data []byte, cli client.TxnClient) (processHelper, error) {
procInfo := &pipeline.ProcessInfo{}
err := procInfo.Unmarshal(data)
if err != nil {
return processHelper{}, err
}
if len(procInfo.GetAnalysisNodeList()) == 0 {
panic(fmt.Sprintf("empty plan: %s", procInfo.Sql))
}
result := processHelper{
id: procInfo.Id,
lim: convertToProcessLimitation(procInfo.Lim),
unixTime: procInfo.UnixTime,
accountId: procInfo.AccountId,
txnClient: cli,
analysisNodeList: procInfo.GetAnalysisNodeList(),
}
result.txnOperator, err = cli.NewWithSnapshot([]byte(procInfo.Snapshot))
if err != nil {
return processHelper{}, err
}
result.sessionInfo, err = convertToProcessSessionInfo(procInfo.SessionInfo)
if err != nil {
return processHelper{}, err
}
return result, nil
}
func (receiver *messageReceiverOnServer) GetProcByUuid(uid uuid.UUID) (*process.Process, error) {
getCtx, getCancel := context.WithTimeout(context.Background(), HandleNotifyTimeout)
var opProc *process.Process
var ok bool
for {
select {
case <-getCtx.Done():
colexec.Srv.GetProcByUuid(uid, true)
getCancel()
return nil, moerr.NewInternalError(receiver.ctx, "get dispatch process by uuid timeout")
case <-receiver.ctx.Done():
colexec.Srv.GetProcByUuid(uid, true)
getCancel()
return nil, nil
default:
if opProc, ok = colexec.Srv.GetProcByUuid(uid, false); !ok {
// it's bad to call the Gosched() here.
// cut the HandleNotifyTimeout to 1ms, 1ms, 2ms, 3ms, 5ms, 8ms..., and use them as waiting time may be a better way.
runtime.Gosched()
} else {
getCancel()
return opProc, nil
}
}
}
}