github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/logstore/driver/logservicedriver/entry.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 "bytes" 19 "encoding/binary" 20 "io" 21 "math" 22 "sync" 23 "sync/atomic" 24 25 "github.com/matrixorigin/matrixone/pkg/common/moerr" 26 "github.com/matrixorigin/matrixone/pkg/logservice" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver" 29 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry" 30 ) 31 32 type MetaType uint8 33 34 const ( 35 TInvalid MetaType = iota 36 TNormal 37 TReplay 38 ) 39 40 type meta struct { 41 metaType MetaType 42 appended uint64 43 addr map[uint64]uint64 44 payloadSize uint64 45 } 46 47 func newMeta() *meta { 48 return &meta{addr: make(map[uint64]uint64), metaType: TNormal} 49 } 50 func (m *meta) SetType(t MetaType) { 51 m.metaType = t 52 } 53 func (m *meta) GetType() MetaType { 54 return m.metaType 55 } 56 func (m *meta) SetAppended(appended uint64) { 57 m.appended = appended 58 } 59 func (m *meta) GetMinLsn() uint64 { 60 min := uint64(0) 61 min = math.MaxUint64 62 for lsn := range m.addr { 63 if lsn < min { 64 min = lsn 65 } 66 } 67 return min 68 } 69 func (m *meta) GetMaxLsn() uint64 { 70 max := uint64(0) 71 for lsn := range m.addr { 72 if lsn > max { 73 max = lsn 74 } 75 } 76 return max 77 } 78 func (m *meta) WriteTo(w io.Writer) (n int64, err error) { 79 if err = binary.Write(w, binary.BigEndian, m.metaType); err != nil { 80 return 81 } 82 n += 1 83 if err = binary.Write(w, binary.BigEndian, m.appended); err != nil { 84 return 85 } 86 n += 8 87 length := uint16(len(m.addr)) 88 if err = binary.Write(w, binary.BigEndian, length); err != nil { 89 return 90 } 91 n += 2 92 for lsn, offset := range m.addr { 93 if err = binary.Write(w, binary.BigEndian, lsn); err != nil { 94 return 95 } 96 n += 8 97 if err = binary.Write(w, binary.BigEndian, offset); err != nil { 98 return 99 } 100 n += 8 101 } 102 if err = binary.Write(w, binary.BigEndian, m.payloadSize); err != nil { 103 return 104 } 105 n += 8 106 return 107 } 108 109 func (m *meta) ReadFrom(r io.Reader) (n int64, err error) { 110 if err = binary.Read(r, binary.BigEndian, &m.metaType); err != nil { 111 return 112 } 113 n += 1 114 if err = binary.Read(r, binary.BigEndian, &m.appended); err != nil { 115 return 116 } 117 n += 8 118 length := uint16(0) 119 if err = binary.Read(r, binary.BigEndian, &length); err != nil { 120 return 121 } 122 n += 2 123 m.addr = make(map[uint64]uint64) 124 for i := 0; i < int(length); i++ { 125 lsn := uint64(0) 126 if err = binary.Read(r, binary.BigEndian, &lsn); err != nil { 127 return 128 } 129 n += 8 130 offset := uint64(0) 131 if err = binary.Read(r, binary.BigEndian, &offset); err != nil { 132 return 133 } 134 n += 8 135 m.addr[lsn] = offset 136 } 137 if err = binary.Read(r, binary.BigEndian, &m.payloadSize); err != nil { 138 return 139 } 140 n += 8 141 return 142 } 143 144 func (m *meta) Unmarshal(buf []byte) error { 145 bbuf := bytes.NewBuffer(buf) 146 _, err := m.ReadFrom(bbuf) 147 return err 148 } 149 150 func (m *meta) Marshal() (buf []byte, err error) { 151 var bbuf bytes.Buffer 152 if _, err = m.WriteTo(&bbuf); err != nil { 153 return 154 } 155 buf = bbuf.Bytes() 156 return 157 } 158 159 // read: logrecord -> meta+payload -> entry 160 // write: entries+meta -> payload -> record 161 type recordEntry struct { 162 *meta 163 entries []*entry.Entry 164 cmd *ReplayCmd 165 166 payload []byte 167 unmarshaled atomic.Uint32 168 mashalMu sync.RWMutex 169 } 170 171 func newRecordEntry() *recordEntry { 172 return &recordEntry{entries: make([]*entry.Entry, 0), meta: newMeta()} 173 } 174 175 func newEmptyRecordEntry(r logservice.LogRecord) *recordEntry { 176 payload := make([]byte, len(r.Payload())) 177 copy(payload, r.Payload()) 178 return &recordEntry{payload: payload, meta: newMeta(), mashalMu: sync.RWMutex{}} 179 } 180 181 func (r *recordEntry) replay(h driver.ApplyHandle) (addr *common.ClosedIntervals) { 182 bbuf := bytes.NewBuffer(r.payload) 183 lsns := make([]uint64, 0) 184 for lsn := range r.meta.addr { 185 lsns = append(lsns, lsn) 186 e := entry.NewEmptyEntry() 187 e.ReadFrom(bbuf) 188 h(e) 189 e.Entry.Free() 190 } 191 intervals := common.NewClosedIntervalsBySlice(lsns) 192 return intervals 193 } 194 func (r *recordEntry) append(e *entry.Entry) { 195 r.entries = append(r.entries, e) 196 r.meta.addr[e.Lsn] = uint64(r.payloadSize) 197 r.payloadSize += uint64(e.GetSize()) 198 } 199 200 func (r *recordEntry) WriteTo(w io.Writer) (n int64, err error) { 201 n1, err := r.meta.WriteTo(w) 202 if err != nil { 203 return 0, err 204 } 205 n += n1 206 switch r.meta.metaType { 207 case TNormal: 208 for _, e := range r.entries { 209 n1, err = e.WriteTo(w) 210 if err != nil { 211 return 212 } 213 n += n1 214 } 215 case TReplay: 216 n1, err = r.cmd.WriteTo(w) 217 if err != nil { 218 return 219 } 220 n += n1 221 default: 222 panic("invalid type") 223 } 224 return 225 } 226 227 func (r *recordEntry) ReadFrom(reader io.Reader) (n int64, err error) { 228 n1, err := r.meta.ReadFrom(reader) 229 if err != nil { 230 return 0, err 231 } 232 n += n1 233 payload := make([]byte, r.meta.payloadSize) 234 n2, err := reader.Read(payload) 235 if err != nil { 236 return 0, err 237 } 238 if n2 != int(r.meta.payloadSize) { 239 panic(moerr.NewInternalErrorNoCtx("logic err: err is %v, expect %d, get %d", err, r.meta.payloadSize, n2)) 240 } 241 r.payload = payload 242 return 243 } 244 245 func (r *recordEntry) Unmarshal(buf []byte) error { 246 bbuf := bytes.NewBuffer(buf) 247 _, err := r.ReadFrom(bbuf) 248 return err 249 } 250 251 func (r *recordEntry) Marshal() (buf []byte, err error) { 252 var bbuf bytes.Buffer 253 if _, err = r.WriteTo(&bbuf); err != nil { 254 return 255 } 256 buf = bbuf.Bytes() 257 return 258 } 259 func (r *recordEntry) prepareRecord() (size int) { 260 var err error 261 r.payload, err = r.Marshal() 262 if err != nil { 263 panic(err) 264 } 265 return len(r.payload) 266 } 267 268 func (r *recordEntry) unmarshal() { 269 if r.unmarshaled.Load() == 1 { 270 return 271 } 272 r.mashalMu.Lock() 273 defer r.mashalMu.Unlock() 274 if r.unmarshaled.Load() == 1 { 275 return 276 } 277 buf := r.payload 278 r.payload = nil 279 err := r.Unmarshal(buf) 280 if err != nil { 281 panic(err) 282 } 283 r.unmarshaled.Store(1) 284 } 285 286 func (r *recordEntry) readEntry(lsn uint64) *entry.Entry { 287 r.unmarshal() 288 offset := r.meta.addr[lsn] 289 bbuf := bytes.NewBuffer(r.payload[offset:]) 290 e := entry.NewEmptyEntry() 291 e.ReadFrom(bbuf) 292 e.Lsn = lsn 293 return e 294 }