github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logstore/driver/logservicedriver/read.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 "sync" 20 "time" 21 22 "github.com/matrixorigin/matrixone/pkg/common/moerr" 23 "github.com/matrixorigin/matrixone/pkg/logservice" 24 "github.com/matrixorigin/matrixone/pkg/logutil" 25 pb "github.com/matrixorigin/matrixone/pkg/pb/logservice" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry" 27 ) 28 29 var ErrRecordNotFound = moerr.NewInternalErrorNoCtx("driver read cache: lsn not found") 30 var ErrAllRecordsRead = moerr.NewInternalErrorNoCtx("driver read cache: all records are read") 31 32 type readCache struct { 33 lsns []uint64 34 records map[uint64]*recordEntry 35 readMu sync.RWMutex 36 } 37 38 func newReadCache() *readCache { 39 return &readCache{ 40 lsns: make([]uint64, 0), 41 records: make(map[uint64]*recordEntry), 42 readMu: sync.RWMutex{}, 43 } 44 } 45 46 func (d *LogServiceDriver) Read(drlsn uint64) (*entry.Entry, error) { 47 lsn, err := d.tryGetLogServiceLsnByDriverLsn(drlsn) 48 if err != nil { 49 panic(err) 50 } 51 d.readMu.RLock() 52 r, err := d.readFromCache(lsn) 53 d.readMu.RUnlock() 54 if err != nil { 55 d.readMu.Lock() 56 r, err = d.readFromCache(lsn) 57 if err == nil { 58 d.readMu.Unlock() 59 return r.readEntry(drlsn), nil 60 } 61 d.readSmallBatchFromLogService(lsn) 62 r, err = d.readFromCache(lsn) 63 if err != nil { 64 logutil.Debugf("try read %d", lsn) 65 panic(err) 66 } 67 d.readMu.Unlock() 68 } 69 return r.readEntry(drlsn), nil 70 } 71 72 func (d *LogServiceDriver) readFromCache(lsn uint64) (*recordEntry, error) { 73 if len(d.records) == 0 { 74 return nil, ErrAllRecordsRead 75 } 76 record, ok := d.records[lsn] 77 if !ok { 78 return nil, ErrRecordNotFound 79 } 80 return record, nil 81 } 82 83 func (d *LogServiceDriver) appendRecords(records []logservice.LogRecord, firstlsn uint64, fn func(uint64, *recordEntry), maxsize int) { 84 lsns := make([]uint64, 0) 85 cnt := 0 86 for i, record := range records { 87 if record.GetType() != pb.UserRecord { 88 continue 89 } 90 lsn := firstlsn + uint64(i) 91 cnt++ 92 if maxsize != 0 { 93 if cnt > maxsize { 94 break 95 } 96 } 97 _, ok := d.records[lsn] 98 if ok { 99 continue 100 } 101 d.records[lsn] = newEmptyRecordEntry(record) 102 lsns = append(lsns, lsn) 103 if fn != nil { 104 fn(lsn, d.records[lsn]) 105 } 106 } 107 d.lsns = append(d.lsns, lsns...) 108 } 109 110 func (d *LogServiceDriver) dropRecords() { 111 drop := len(d.lsns) - d.config.ReadCacheSize 112 lsns := d.lsns[:drop] 113 for _, lsn := range lsns { 114 delete(d.records, lsn) 115 } 116 d.lsns = d.lsns[drop:] 117 } 118 func (d *LogServiceDriver) dropRecordByLsn(lsn uint64) { 119 delete(d.records, lsn) 120 } 121 122 func (d *LogServiceDriver) resetReadCache() { 123 for lsn := range d.records { 124 delete(d.records, lsn) 125 } 126 } 127 128 func (d *LogServiceDriver) readSmallBatchFromLogService(lsn uint64) { 129 _, records := d.readFromLogService(lsn, int(d.config.RecordSize)) 130 if len(records) == 0 { 131 _, records = d.readFromLogService(lsn, MaxReadSize) 132 } 133 d.appendRecords(records, lsn, nil, 1) 134 if !d.IsReplaying() && len(d.lsns) > d.config.ReadCacheSize { 135 d.dropRecords() 136 } 137 } 138 139 func (d *LogServiceDriver) readFromLogServiceInReplay(lsn uint64, size int, fn func(lsn uint64, r *recordEntry)) (uint64, uint64) { 140 nextLsn, records := d.readFromLogService(lsn, size) 141 safeLsn := uint64(0) 142 d.appendRecords(records, lsn, func(lsn uint64, r *recordEntry) { 143 r.unmarshal() 144 if safeLsn < r.appended { 145 safeLsn = r.appended 146 } 147 fn(lsn, r) 148 }, 0) 149 return nextLsn, safeLsn 150 } 151 152 func (d *LogServiceDriver) readFromLogService(lsn uint64, size int) (nextLsn uint64, records []logservice.LogRecord) { 153 client, err := d.clientPool.Get() 154 defer d.clientPool.Put(client) 155 if err != nil { 156 panic(err) 157 } 158 t0 := time.Now() 159 ctx, cancel := context.WithTimeout(context.Background(), d.config.ReadDuration) 160 records, nextLsn, err = client.c.Read(ctx, lsn, uint64(size)) 161 cancel() 162 if err != nil { 163 err = RetryWithTimeout(d.config.RetryTimeout, func() (shouldReturn bool) { 164 logutil.Infof("LogService Driver: retry read err is %v", err) 165 ctx, cancel := context.WithTimeout(context.Background(), d.config.ReadDuration) 166 records, nextLsn, err = client.c.Read(ctx, lsn, uint64(size)) 167 cancel() 168 return err == nil 169 }) 170 if err != nil { 171 panic(err) 172 } 173 } 174 d.readDuration += time.Since(t0) 175 return 176 }