github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logstore/driver/logservicedriver/appender.go (about) 1 // Copyright 2021 Matrix Origin 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package logservicedriver 16 17 import ( 18 "context" 19 gotrace "runtime/trace" 20 "sync" 21 "time" 22 23 "github.com/matrixorigin/matrixone/pkg/logutil" 24 v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 25 "github.com/matrixorigin/matrixone/pkg/util/trace" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry" 27 ) 28 29 type driverAppender struct { 30 client *clientWithRecord 31 appendlsn uint64 32 logserviceLsn uint64 33 entry *recordEntry 34 contextDuration time.Duration 35 wg sync.WaitGroup //wait client 36 } 37 38 func newDriverAppender() *driverAppender { 39 return &driverAppender{ 40 entry: newRecordEntry(), 41 wg: sync.WaitGroup{}, 42 } 43 } 44 45 func (a *driverAppender) appendEntry(e *entry.Entry) { 46 a.entry.append(e) 47 } 48 49 func (a *driverAppender) append(retryTimout, appendTimeout time.Duration) { 50 _, task := gotrace.NewTask(context.Background(), "logservice.append") 51 start := time.Now() 52 defer func() { 53 v2.LogTailAppendDurationHistogram.Observe(time.Since(start).Seconds()) 54 task.End() 55 }() 56 57 size := a.entry.prepareRecord() 58 // if size > int(common.K)*20 { //todo 59 // panic(moerr.NewInternalError("record size %d, larger than max size 20K", size)) 60 // } 61 a.client.TryResize(size) 62 logutil.Debugf("Log Service Driver: append start prepare %p", a.client.record.Data) 63 record := a.client.record 64 copy(record.Payload(), a.entry.payload) 65 record.ResizePayload(size) 66 defer logSlowAppend()() 67 ctx, cancel := context.WithTimeout(context.Background(), appendTimeout) 68 69 var timeoutSpan trace.Span 70 // Before issue#10467 is resolved, we skip this span, 71 // avoiding creating too many goroutines, which affects the performance. 72 ctx, timeoutSpan = trace.Debug(ctx, "appender", 73 trace.WithProfileGoroutine(), 74 trace.WithProfileHeap(), 75 trace.WithProfileCpuSecs(time.Second*10)) 76 defer timeoutSpan.End() 77 78 v2.LogTailBytesHistogram.Observe(float64(size)) 79 logutil.Debugf("Log Service Driver: append start %p", a.client.record.Data) 80 lsn, err := a.client.c.Append(ctx, record) 81 if err != nil { 82 logutil.Errorf("append failed: %v", err) 83 } 84 cancel() 85 if err != nil { 86 err = RetryWithTimeout(retryTimout, func() (shouldReturn bool) { 87 ctx, cancel := context.WithTimeout(context.Background(), appendTimeout) 88 ctx, timeoutSpan = trace.Debug(ctx, "appender retry", 89 trace.WithProfileGoroutine(), 90 trace.WithProfileHeap(), 91 trace.WithProfileCpuSecs(time.Second*10)) 92 defer timeoutSpan.End() 93 lsn, err = a.client.c.Append(ctx, record) 94 cancel() 95 if err != nil { 96 logutil.Errorf("append failed: %v", err) 97 } 98 return err == nil 99 }) 100 } 101 logutil.Debugf("Log Service Driver: append end %p", a.client.record.Data) 102 if err != nil { 103 logutil.Infof("size is %d", size) 104 logutil.Panic(err.Error()) 105 } 106 a.logserviceLsn = lsn 107 a.wg.Done() 108 } 109 110 func (a *driverAppender) waitDone() { 111 a.wg.Wait() 112 } 113 114 func (a *driverAppender) freeEntries() { 115 for _, e := range a.entry.entries { 116 e.DoneWithErr(nil) 117 } 118 } 119 120 func logSlowAppend() func() { 121 const slowAppend = 1 * time.Second 122 start := time.Now() 123 return func() { 124 elapsed := time.Since(start) 125 if elapsed >= slowAppend { 126 logutil.Warnf("append to logservice took %s", elapsed) 127 } 128 } 129 }