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