github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logstore/entry/base.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 entry 16 17 import ( 18 "bytes" 19 "fmt" 20 "io" 21 "os" 22 "sync" 23 "time" 24 25 "github.com/matrixorigin/matrixone/pkg/container/types" 26 "github.com/matrixorigin/matrixone/pkg/logutil" 27 "github.com/matrixorigin/matrixone/pkg/objectio" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 29 ) 30 31 var ( 32 _basePool = sync.Pool{New: func() any { 33 return &Base{ 34 descriptor: newDescriptor(), 35 } 36 }} 37 ) 38 39 type CkpRanges struct { 40 Group uint32 41 Ranges *common.ClosedIntervals 42 Command map[uint64]CommandInfo 43 } 44 45 func (r CkpRanges) String() string { 46 s := fmt.Sprintf("G%d-%v", r.Group, r.Ranges) 47 for lsn, cmd := range r.Command { 48 s = fmt.Sprintf("%s[%d-%v/%d]", s, lsn, cmd.CommandIds, cmd.Size) 49 } 50 return s 51 } 52 53 type CommandInfo struct { 54 CommandIds []uint32 55 Size uint32 56 } 57 type Info struct { 58 Group uint32 59 Checkpoints []*CkpRanges 60 GroupLSN uint64 61 62 TargetLsn uint64 63 Info any 64 } 65 66 func NewEmptyInfo() *Info { 67 return &Info{} 68 } 69 func (info *Info) WriteTo(w io.Writer) (n int64, err error) { 70 if _, err = w.Write(types.EncodeUint32(&info.Group)); err != nil { 71 return 72 } 73 n += 4 74 if _, err = w.Write(types.EncodeUint64(&info.GroupLSN)); err != nil { 75 return 76 } 77 n += 8 78 if _, err = w.Write(types.EncodeUint64(&info.TargetLsn)); err != nil { 79 return 80 } 81 n += 8 82 length := uint64(len(info.Checkpoints)) 83 if _, err = w.Write(types.EncodeUint64(&length)); err != nil { 84 return 85 } 86 n += 8 87 for _, ckps := range info.Checkpoints { 88 if _, err = w.Write(types.EncodeUint32(&ckps.Group)); err != nil { 89 return 90 } 91 n += 4 92 var n2 int64 93 n2, err = ckps.Ranges.WriteTo(w) 94 if err != nil { 95 return 96 } 97 n += n2 98 cmdLength := uint64(len(ckps.Command)) 99 if _, err = w.Write(types.EncodeUint64(&cmdLength)); err != nil { 100 return 101 } 102 n += 8 103 for lsn, cmd := range ckps.Command { 104 if _, err = w.Write(types.EncodeUint64(&lsn)); err != nil { 105 return 106 } 107 n += 8 108 cmdIdxLength := uint32(len(cmd.CommandIds)) 109 if _, err = w.Write(types.EncodeUint32(&cmdIdxLength)); err != nil { 110 return 111 } 112 n += 4 113 for _, id := range cmd.CommandIds { 114 if _, err = w.Write(types.EncodeUint32(&id)); err != nil { 115 return 116 } 117 n += 4 118 } 119 if _, err = w.Write(types.EncodeUint32(&cmd.Size)); err != nil { 120 return 121 } 122 n += 4 123 } 124 } 125 return 126 } 127 func (info *Info) Marshal() (buf []byte, err error) { 128 var bbuf bytes.Buffer 129 if _, err = info.WriteTo(&bbuf); err != nil { 130 return 131 } 132 buf = bbuf.Bytes() 133 return 134 } 135 func (info *Info) ReadFrom(r io.Reader) (n int64, err error) { 136 if _, err = r.Read(types.EncodeUint32(&info.Group)); err != nil { 137 return 138 } 139 n += 4 140 if _, err = r.Read(types.EncodeUint64(&info.GroupLSN)); err != nil { 141 return 142 } 143 n += 8 144 if _, err = r.Read(types.EncodeUint64(&info.TargetLsn)); err != nil { 145 return 146 } 147 n += 8 148 length := uint64(0) 149 if _, err = r.Read(types.EncodeUint64(&length)); err != nil { 150 return 151 } 152 n += 8 153 info.Checkpoints = make([]*CkpRanges, length) 154 for i := 0; i < int(length); i++ { 155 ckps := &CkpRanges{} 156 if _, err = r.Read(types.EncodeUint32(&ckps.Group)); err != nil { 157 return 158 } 159 n += 4 160 ckps.Ranges = common.NewClosedIntervals() 161 var n2 int64 162 n2, err = ckps.Ranges.ReadFrom(r) 163 if err != nil { 164 return 165 } 166 n += n2 167 cmdLength := uint64(0) 168 if _, err = r.Read(types.EncodeUint64(&cmdLength)); err != nil { 169 return 170 } 171 n += 8 172 ckps.Command = make(map[uint64]CommandInfo) 173 for i := 0; i < int(cmdLength); i++ { 174 lsn := uint64(0) 175 if _, err = r.Read(types.EncodeUint64(&lsn)); err != nil { 176 return 177 } 178 n += 8 179 cmd := &CommandInfo{} 180 cmdIdxLength := uint32(0) 181 if _, err = r.Read(types.EncodeUint32(&cmdIdxLength)); err != nil { 182 return 183 } 184 n += 4 185 cmd.CommandIds = make([]uint32, cmdIdxLength) 186 for i := 0; i < int(cmdIdxLength); i++ { 187 if _, err = r.Read(types.EncodeUint32(&cmd.CommandIds[i])); err != nil { 188 return 189 } 190 n += 4 191 } 192 if _, err = r.Read(types.EncodeUint32(&cmd.Size)); err != nil { 193 return 194 } 195 n += 4 196 ckps.Command[lsn] = *cmd 197 } 198 info.Checkpoints[i] = ckps 199 } 200 return 201 } 202 func (info *Info) Unmarshal(buf []byte) error { 203 bbuf := bytes.NewBuffer(buf) 204 _, err := info.ReadFrom(bbuf) 205 return err 206 } 207 208 func (info *Info) ToString() string { 209 switch info.Group { 210 case GTCKp: 211 s := "checkpoint entry" 212 for _, ranges := range info.Checkpoints { 213 s = fmt.Sprintf("%s%s", s, ranges) 214 } 215 s = fmt.Sprintf("%s\n", s) 216 return s 217 default: 218 s := fmt.Sprintf("customized entry G%d<%d>", info.Group, info.GroupLSN) 219 s = fmt.Sprintf("%s\n", s) 220 return s 221 } 222 } 223 224 type Base struct { 225 *descriptor 226 payload []byte 227 info any 228 infobuf []byte 229 wg sync.WaitGroup 230 t0 time.Time 231 printTime bool 232 err error 233 } 234 235 func GetBase() *Base { 236 b := _basePool.Get().(*Base) 237 if b.GetPayloadSize() != 0 { 238 logutil.Debugf("payload size is %d", b.GetPayloadSize()) 239 panic("wrong payload size") 240 } 241 b.wg.Add(1) 242 return b 243 } 244 func (b *Base) StartTime() { 245 b.t0 = time.Now() 246 } 247 func (b *Base) Duration() time.Duration { 248 return time.Since(b.t0) 249 } 250 func (b *Base) PrintTime() { 251 b.printTime = true 252 } 253 func (b *Base) IsPrintTime() bool { 254 return b.printTime 255 } 256 func (b *Base) reset() { 257 b.descriptor.reset() 258 b.payload = nil 259 b.info = nil 260 b.infobuf = nil 261 b.wg = sync.WaitGroup{} 262 b.t0 = time.Time{} 263 b.printTime = false 264 b.err = nil 265 } 266 func (b *Base) GetInfoBuf() []byte { 267 return b.infobuf 268 } 269 func (b *Base) SetInfoBuf(buf []byte) { 270 b.SetInfoSize(len(buf)) 271 b.infobuf = buf 272 } 273 func (b *Base) GetError() error { 274 return b.err 275 } 276 277 func (b *Base) WaitDone() error { 278 b.wg.Wait() 279 return b.err 280 } 281 282 func (b *Base) DoneWithErr(err error) { 283 b.err = err 284 b.wg.Done() 285 } 286 287 func (b *Base) Free() { 288 b.reset() 289 if b.GetPayloadSize() != 0 { 290 logutil.Debugf("payload size is %d", b.GetPayloadSize()) 291 panic("wrong payload size") 292 } 293 _basePool.Put(b) 294 } 295 296 func (b *Base) GetPayload() []byte { 297 return b.payload 298 } 299 300 func (b *Base) SetInfo(info any) { 301 b.info = info 302 } 303 304 func (b *Base) GetInfo() any { 305 return b.info 306 } 307 308 func (b *Base) UnmarshalFromNode(n []byte, own bool) error { 309 if own { 310 b.payload = n 311 } else { 312 copy(b.payload, n) 313 } 314 b.SetPayloadSize(len(b.payload)) 315 return nil 316 } 317 318 func (b *Base) SetPayload(buf []byte) error { 319 b.payload = buf 320 b.SetPayloadSize(len(buf)) 321 return nil 322 } 323 324 func (b *Base) Unmarshal(buf []byte) error { 325 bbuf := bytes.NewBuffer(buf) 326 _, err := b.ReadFrom(bbuf) 327 return err 328 } 329 func (b *Base) GetLsn() (gid uint32, lsn uint64) { 330 v := b.GetInfo() 331 if v == nil { 332 return 333 } 334 info := v.(*Info) 335 gid = info.Group 336 lsn = info.GroupLSN 337 return 338 } 339 func (b *Base) Marshal() (buf []byte, err error) { 340 var bbuf bytes.Buffer 341 if _, err = b.WriteTo(&bbuf); err != nil { 342 return 343 } 344 buf = bbuf.Bytes() 345 return 346 } 347 348 func (b *Base) ReadFrom(r io.Reader) (int64, error) { 349 metaBuf := b.GetMetaBuf() 350 _, err := r.Read(metaBuf) 351 if err != nil { 352 return 0, err 353 } 354 355 b.payload = make([]byte, b.GetPayloadSize()) 356 n1 := 0 357 if b.GetInfoSize() != 0 { 358 infoBuf := make([]byte, b.GetInfoSize()) 359 n, err := r.Read(infoBuf) 360 n1 += n 361 if err != nil { 362 return int64(n1), err 363 } 364 head := objectio.DecodeIOEntryHeader(b.descBuf) 365 codec := objectio.GetIOEntryCodec(*head) 366 vinfo, err := codec.Decode(infoBuf) 367 info := vinfo.(*Info) 368 if err != nil { 369 return int64(n1), err 370 } 371 b.SetInfo(info) 372 } 373 n2, err := r.Read(b.payload) 374 if err != nil { 375 return int64(n2), err 376 } 377 return int64(n1 + n2), nil 378 } 379 380 func (b *Base) UnmarshalBinary(buf []byte) (n int64, err error) { 381 copy(b.GetMetaBuf(), buf[:b.GetMetaSize()]) 382 n += int64(b.GetMetaSize()) 383 384 infoSize := b.GetInfoSize() 385 if infoSize != 0 { 386 head := objectio.DecodeIOEntryHeader(b.descBuf) 387 codec := objectio.GetIOEntryCodec(*head) 388 vinfo, err := codec.Decode(buf[n : n+int64(infoSize)]) 389 info := vinfo.(*Info) 390 if err != nil { 391 return n, err 392 } 393 b.SetInfo(info) 394 n += int64(infoSize) 395 } 396 397 payloadSize := b.GetPayloadSize() 398 b.payload = buf[n : n+int64(payloadSize)] 399 n += int64(payloadSize) 400 return 401 } 402 403 func (b *Base) ReadAt(r *os.File, offset int) (int, error) { 404 metaBuf := b.GetMetaBuf() 405 n, err := r.ReadAt(metaBuf, int64(offset)) 406 if err != nil { 407 return n, err 408 } 409 b.payload = make([]byte, b.GetPayloadSize()) 410 411 offset += len(b.GetMetaBuf()) 412 infoBuf := make([]byte, b.GetInfoSize()) 413 n1, err := r.ReadAt(infoBuf, int64(offset)) 414 if err != nil { 415 return n + n1, err 416 } 417 offset += n1 418 b.SetInfoBuf(infoBuf) 419 420 head := objectio.DecodeIOEntryHeader(b.descBuf) 421 codec := objectio.GetIOEntryCodec(*head) 422 vinfo, err := codec.Decode(infoBuf) 423 info := vinfo.(*Info) 424 if err != nil { 425 return n + n1, err 426 } 427 b.SetInfo(info) 428 n2, err := r.ReadAt(b.payload, int64(offset)) 429 if err != nil { 430 return n2, err 431 } 432 return n + n1 + n2, nil 433 } 434 435 func (b *Base) PrepareWrite() { 436 if b.info == nil { 437 return 438 } 439 buf, err := b.info.(*Info).Marshal() 440 if err != nil { 441 panic(err) 442 } 443 b.SetInfoBuf(buf) 444 } 445 446 func (b *Base) WriteTo(w io.Writer) (int64, error) { 447 b.descriptor.SetVersion(IOET_WALEntry_CurrVer) 448 n1, err := b.descriptor.WriteTo(w) 449 if err != nil { 450 return n1, err 451 } 452 n2, err := w.Write(b.GetInfoBuf()) 453 if err != nil { 454 return int64(n2), err 455 } 456 n3, err := w.Write(b.payload) 457 if err != nil { 458 return int64(n3), err 459 } 460 return n1 + int64(n2) + int64(n3), err 461 }